NIFI-15483: Route PublishAMQP to failure on undeliverable messages instead of silent success#11213
Conversation
5da2156 to
ace38df
Compare
|
@rakesh-rsky Thanks for working on this issue. Please note the unit tests are failing. I tried the new error handling in my local environment but I'm getting the following runtime error (instead of routing the FlowFile to failure): |
|
@turcsanyip Thank you for validating this. Root cause: When the exchange does not exist, the broker closes the channel with This has been fixed in the latest commits — |
46fe897 to
89f5fe6
Compare
…r cannot deliver message PublishAMQP uses mandatory=true on basicPublish() so the broker returns messages it cannot route to any queue. However, the return arrives asynchronously via ReturnListener.handleReturn() on the AMQP I/O thread while the publishing thread had already moved on to session.transfer(REL_SUCCESS). The UndeliverableMessageLogger only logged a warning — it never signaled failure back to publish() or onTrigger(), so every unroutable message was silently counted as a success despite never reaching any consumer. Fix: - Enabled Publisher Confirms (channel.confirmSelect()) in the constructor. The broker's basic.return frame for an unroutable message is guaranteed to arrive before the corresponding confirm frame, so waitForConfirms() acts as a synchronization barrier that makes return detection reliable. - Added an AtomicReference<String> field (undeliverableReturnReason) that UndeliverableMessageLogger.handleReturn() populates with exchange/routingKey/ replyCode/replyText when a message is returned. - publish() now: resets the field before each call, calls waitForConfirms(5s) to synchronize with the broker, then checks the field and throws AMQPException if the message was returned — causing onTrigger() to route to REL_FAILURE. - Broker NACKs (e.g., resource alarm) are also now surfaced as AMQPException because waitForConfirms() returns false on NACK. - Added regression tests to verify that AMQPPublisher and PublishAMQP correctly route FlowFiles to REL_FAILURE for all broker-side failure modes: - Added ShutdownSignalException to the catch block in AMQPPublisher.publish() - Converts the channel-close signal into AMQPException so PublishAMQP routes the FlowFile to REL_FAILURE with a descriptive error message - Added ShutdownSignalException import Co-authored-by: Rakesh Kumar Singh <rsky.rakesh@gmail.com>
89f5fe6 to
99cf2e5
Compare
|
@rakesh-rsky I checked the latest changes and the error handling is correct now functionally. However, the synchronous wait led to significant performance degradation compared to the original version. According to my measurement:
So even if the error handling is improved, I don't think we can add the change as-is due to the performance effect. I suggest adding a feature flag property (e.g. |
Summary
PublishAMQPsilently routes FlowFiles toREL_SUCCESSeven when the AMQP broker cannot deliver the message, causing silent data loss.Two failure modes are addressed:
Undeliverable message (
basic.return) — broker returns the message when no queue is bound to the exchange/routing-key. The fix uses AMQP Publisher Confirms +basic.returnto detect and surface delivery failures, routing the FlowFile toREL_FAILURE.Exchange not found (
ShutdownSignalException) — when the exchange does not exist, the broker closes the channel with404 NOT_FOUND, causingwaitForConfirms()to throwShutdownSignalException. This is now caught and converted toAMQPExceptionso the FlowFile routes toREL_FAILUREinstead of causing an unhandled processor failure.Testing
failurewith the broker's return reasonfailureinstead of unhandled processor errorsuccessFixes: https://issues.apache.org/jira/browse/NIFI-15483